0be36b
@@ -19,6 +19,7 @@
 
 import java.io.IOException;
 import java.net.URI;
+import java.util.Iterator;
 import java.util.List;
 
 import org.apache.commons.lang.StringUtils;
@@ -27,6 +28,7 @@
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.InvalidObjectException;
 import org.apache.hadoop.hive.metastore.api.InvalidOperationException;
 import org.apache.hadoop.hive.metastore.api.MetaException;
@@ -97,10 +99,14 @@
public void alterTable(RawStore msdb, Warehouse wh, String dbname,
             + newt.getTableName() + " doesn't exist");
       }
 
-      // check that partition keys have not changed, except for virtual views
+      //check that partition keys have not changed, except for virtual views
+      //however, allow the partition comments to change
+      boolean partKeysPartiallyEqual = checkPartialPartKeysEqual(oldt.getPartitionKeys(),
+          newt.getPartitionKeys());
+
       if(!oldt.getTableType().equals(TableType.VIRTUAL_VIEW.toString())){
         if (oldt.getPartitionKeys().size() != newt.getPartitionKeys().size()
-            || !oldt.getPartitionKeys().containsAll(newt.getPartitionKeys())) {
+            || !partKeysPartiallyEqual) {
           throw new InvalidOperationException(
               "partition keys can not be changed.");
         }
@@ -215,4 +221,29 @@
public void alterTable(RawStore msdb, Warehouse wh, String dbname,
       }
     }
   }
-}
+
+  private boolean checkPartialPartKeysEqual(List<FieldSchema> oldPartKeys,
+      List<FieldSchema> newPartKeys) {
+    //return true if both are null, or false if one is null and the other isn't
+    if (newPartKeys == null || oldPartKeys == null) {
+      return oldPartKeys == newPartKeys;
+    }
+    if (oldPartKeys.size() != newPartKeys.size()) {
+      return false;
+    }
+    Iterator<FieldSchema> oldPartKeysIter = oldPartKeys.iterator();
+    Iterator<FieldSchema> newPartKeysIter = newPartKeys.iterator();
+    FieldSchema oldFs;
+    FieldSchema newFs;
+    while (oldPartKeysIter.hasNext()) {
+      oldFs = oldPartKeysIter.next();
+      newFs = newPartKeysIter.next();
+      if (!oldFs.getName().equals(newFs.getName()) ||
+          !oldFs.getType().equals(newFs.getType())) {
+        return false;
+      }
+    }
+
+    return true;
+  }
+}
\ No newline at end of file
